637224
@@ -61,8 +61,10 @@
 import org.apache.hadoop.fs.FileChecksum;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hive.common.FileUtils;
 import org.apache.hadoop.hive.common.HiveStatsUtils;
 import org.apache.hadoop.hive.common.JavaUtils;
@@ -3670,7 +3672,15 @@
public static boolean moveFile(final HiveConf conf, Path srcf, final Path destf,
                   "Unable to move source " + srcStatus.getPath() + " to destination " + destFile;
 
               if (null == pool) {
-                if(!destFs.rename(srcStatus.getPath(), destFile)) {
+                boolean success = false;
+                if (destFs instanceof DistributedFileSystem) {
+                  ((DistributedFileSystem)destFs).rename(srcStatus.getPath(), destFile, Options.Rename.OVERWRITE);
+                  success = true;
+                } else {
+                  destFs.delete(destFile, false);
+                  success = destFs.rename(srcStatus.getPath(), destFile);
+                }
+                if(!success) {
                   throw new IOException("rename for src path: " + srcStatus.getPath() + " to dest:"
                       + destf + " returned false");
                 }
@@ -3681,7 +3691,15 @@
public Void call() throws HiveException {
                     SessionState.setCurrentSessionState(parentSession);
                     final String group = srcStatus.getGroup();
                     try {
-                      if (!destFs.rename(srcStatus.getPath(), destFile)) {
+                      boolean success = false;
+                      if (destFs instanceof DistributedFileSystem) {
+                        ((DistributedFileSystem)destFs).rename(srcStatus.getPath(), destFile, Options.Rename.OVERWRITE);
+                        success = true;
+                      } else {
+                        destFs.delete(destFile, false);
+                        success = destFs.rename(srcStatus.getPath(), destFile);
+                      }
+                      if (!success) {
                         throw new IOException(
                             "rename for src path: " + srcStatus.getPath() + " to dest path:"
                                 + destFile + " returned false");
